NIFI-14472: Fix potential NPE in PutKinesisFirehose batch accumulation#11211
Conversation
36af390 to
67c2d4c
Compare
|
@rakesh-rsky It looks like the last update to this branch brought in a commit instead of rebasing, can you rebase your branch against |
exceptionfactory
left a comment
There was a problem hiding this comment.
Thanks for addressing this problem @rakesh-rsky. The core functional change appears to be correct, but the comments are unnecessary and the implementation line should be broken out into several lines for readability. If you can make that change, remove the extra commit, and rebase against the current main branch, this should be good to go.
| // Use a single computeIfAbsent().add() call so the list lookup is atomic. | ||
| // The previous two-step pattern (computeIfAbsent then a separate get()) could | ||
| // throw NullPointerException when firehoseStreamName evaluates to null via | ||
| // expression language, because get(null) returned null after the absent-key | ||
| // entry was never actually inserted. (NIFI-14472) | ||
| session.read(flowFile, in -> recordHash.computeIfAbsent(firehoseStreamName, k -> new ArrayList<>()) | ||
| .add(Record.builder().data(SdkBytes.fromInputStream(in)).build())); |
There was a problem hiding this comment.
Commenting on the previous behavior is unnecessary and should be removed. In addition, it would be helpful to separate the callback implementation across multiple lines for readability.
0f4ced8 to
8bf55d0
Compare
exceptionfactory
left a comment
There was a problem hiding this comment.
Thanks for the update @rakesh-rsky, the functional change looks good, but the branch still includes the unrelated change to KubernetesConfigMapStateProviderTest.
8bf55d0 to
f3a3b19
Compare
…eProvider teardown - Collapsed two-step computeIfAbsent/get into a single chained call in PutKinesisFirehose to eliminate the redundant map lookup. - Guarded KubernetesConfigMapStateProvider.shutdown() against null kubernetesClient and logger so it is safe to call on an uninitialized provider.
f3a3b19 to
d9cf185
Compare
exceptionfactory
left a comment
There was a problem hiding this comment.
Thanks again for making the changes @rakesh-rsky, the latest version looks good. +1 merging
Summary
Fixes a race condition / NPE in PutKinesisFirehose.onTrigger() where a two-step computeIfAbsent() + get() on the batch-accumulation map could return null if the entry was evicted between the two calls.
Changes
Testing
Fixes: https://issues.apache.org/jira/browse/NIFI-14472